f8dd99
@@ -521,6 +521,7 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
     final FlushResult result; // indicating a failure result from prepare
     final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
     final TreeMap<byte[], List<Path>> committedFiles;
+    final TreeMap<byte[], Long> storeFlushableSize;
     final long startTime;
     final long flushOpSeqId;
     final long flushedSeqId;
@@ -528,26 +529,29 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
 
     /** Constructs an early exit case */
     PrepareFlushResult(FlushResult result, long flushSeqId) {
-      this(result, null, null, Math.max(0, flushSeqId), 0, 0, 0);
+      this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, 0);
     }
 
     /** Constructs a successful prepare flush result */
     PrepareFlushResult(
       TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
-      TreeMap<byte[], List<Path>> committedFiles, long startTime, long flushSeqId,
+      TreeMap<byte[], List<Path>> committedFiles,
+      TreeMap<byte[], Long> storeFlushableSize, long startTime, long flushSeqId,
       long flushedSeqId, long totalFlushableSize) {
-      this(null, storeFlushCtxs, committedFiles, startTime,
+      this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
         flushSeqId, flushedSeqId, totalFlushableSize);
     }
 
     private PrepareFlushResult(
       FlushResult result,
       TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
-      TreeMap<byte[], List<Path>> committedFiles, long startTime, long flushSeqId,
+      TreeMap<byte[], List<Path>> committedFiles,
+      TreeMap<byte[], Long> storeFlushableSize, long startTime, long flushSeqId,
       long flushedSeqId, long totalFlushableSize) {
       this.result = result;
       this.storeFlushCtxs = storeFlushCtxs;
       this.committedFiles = committedFiles;
+      this.storeFlushableSize = storeFlushableSize;
       this.startTime = startTime;
       this.flushOpSeqId = flushSeqId;
       this.flushedSeqId = flushedSeqId;
@@ -2156,6 +2160,8 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
       = new TreeMap<byte[], StoreFlushContext>(Bytes.BYTES_COMPARATOR);
     TreeMap<byte[], List<Path>> committedFiles = new TreeMap<byte[], List<Path>>(
         Bytes.BYTES_COMPARATOR);
+    TreeMap<byte[], Long> storeFlushableSize
+        = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
     // The sequence id of this flush operation which is used to log FlushMarker and pass to
     // createFlushContext to use as the store file's sequence id. It can be in advance of edits
     // still in the memstore, edits that are in other column families yet to be flushed.
@@ -2194,6 +2200,7 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
           totalFlushableSizeOfFlushableStores += s.getFlushableSize();
           storeFlushCtxs.put(s.getFamily().getName(), s.createFlushContext(flushOpSeqId));
           committedFiles.put(s.getFamily().getName(), null); // for writing stores to WAL
+          storeFlushableSize.put(s.getFamily().getName(), s.getFlushableSize());
         }
 
         // write the snapshot start to WAL
@@ -2260,7 +2267,7 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         mvcc.advanceMemstore(w);
       }
     }
-    return new PrepareFlushResult(storeFlushCtxs, committedFiles, startTime, flushOpSeqId,
+    return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime, flushOpSeqId,
       flushedSeqId, totalFlushableSizeOfFlushableStores);
   }
 
@@ -2335,7 +2342,13 @@
public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
         if (needsCompaction) {
           compactionRequested = true;
         }
-        committedFiles.put(it.next().getFamily().getName(), flush.getCommittedFiles());
+        byte[] storeName = it.next().getFamily().getName();
+        List<Path> storeCommittedFiles = flush.getCommittedFiles();
+        committedFiles.put(storeName, storeCommittedFiles);
+        // Flush committed no files, indicating flush is empty or flush was canceled
+        if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) {
+          totalFlushableSizeOfFlushableStores -= prepareResult.storeFlushableSize.get(storeName);
+        }
       }
       storeFlushCtxs.clear();
 
